839dec
@@ -21,7 +21,6 @@
package org.apache.hadoop.hbase.master;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -40,7 +39,6 @@
import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
-import com.google.common.collect.LinkedHashMultimap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -85,6 +83,8 @@
import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.KeeperException.NodeExistsException;
 import org.apache.zookeeper.data.Stat;
 
+import com.google.common.collect.LinkedHashMultimap;
+
 /**
  * Manages and performs region assignment.
  * <p>
@@ -1061,13 +1061,27 @@
public class AssignmentManager extends ZooKeeperListener {
               ZKUtil.listChildrenAndWatchForNewChildren(
                 watcher, watcher.assignmentZNode);
             if (children != null) {
+              Stat stat = new Stat();
               for (String child : children) {
                 // if region is in transition, we already have a watch
                 // on it, so no need to watch it again. So, as I know for now,
                 // this is needed to watch splitting nodes only.
                 if (!regionStates.isRegionInTransition(child)) {
-                  ZKUtil.watchAndCheckExists(watcher,
-                    ZKUtil.joinZNode(watcher.assignmentZNode, child));
+                  stat.setVersion(0);
+                  byte[] data = ZKAssign.getDataAndWatch(watcher,
+                    ZKUtil.joinZNode(watcher.assignmentZNode, child), stat);
+                  if (data != null && stat.getVersion() > 0) {
+                    try {
+                      RegionTransition rt = RegionTransition.parseFrom(data);
+
+                      //See HBASE-7551, handle splitting too, in case we miss the node change event
+                      if (rt.getEventType() == EventType.RS_ZK_REGION_SPLITTING) {
+                        handleRegion(rt, stat.getVersion());
+                      }
+                    } catch (DeserializationException de) {
+                      LOG.error("error getting data for " + child, de);
+                    }
+                  }
                 }
               }
             }
